Skip to content

[Data] Store _source_paths in object store rather than self to prevent excessive spilling during read task serialization#59999

Merged
bveeramani merged 4 commits intoray-project:masterfrom
pinterest:jiunyulee/revert
Feb 3, 2026
Merged

[Data] Store _source_paths in object store rather than self to prevent excessive spilling during read task serialization#59999
bveeramani merged 4 commits intoray-project:masterfrom
pinterest:jiunyulee/revert

Conversation

@lee1258561
Copy link
Contributor

@lee1258561 lee1258561 commented Jan 9, 2026

Description

This PR reintroducing the issue that A previous PR trying to solve: #55978

Specifically, this add a full list of paths to self and for Datasource self is captured every read_task_fn during serialization and causing this data being duplicated and cause excessive spilling.

We face the similar warning and have spilling behavior:

The serialized size of your read function named 'read_task_fn' is 9.7MB. This size relatively large. As a result, Ray might excessively spill objects during execution. To fix this issue, avoid accessing `self` or other large objects in 'read_task_fn'.

Revert can solve the issue but this attribute seems to have other dependancy.

This PR move the _source_paths to object store during init time and only keep the object ref in self. and introduce a property function to materialize raw path data when needed

Related issues

#55978

Additional information

Optional: Add implementation details, API changes, usage examples, screenshots, etc.

@lee1258561 lee1258561 requested a review from a team as a code owner January 9, 2026 08:30
Copy link
Contributor

@gemini-code-assist gemini-code-assist bot left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Code Review

This pull request reverts a change that added _source_paths to FileBasedDatasource and ParquetDatasource for lineage tracking. As you've noted in the description, this change introduced a significant performance regression due to the increased serialization size of read tasks, leading to excessive object spilling. The revert correctly removes the problematic _source_paths attribute and the corresponding test assertions. This is a necessary fix for the performance issue, and I approve of this change. Given that this re-introduces the issue with lineage tracking, it would be beneficial to create a follow-up ticket to explore a more performant solution for tracking source paths.

@ray-gardener ray-gardener bot added data Ray Data-related issues community-contribution Contributed by the community labels Jan 9, 2026
@lee1258561 lee1258561 added the go add ONLY when ready to merge, run all tests label Jan 9, 2026
Copy link
Contributor

@goutamvenkat-anyscale goutamvenkat-anyscale left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for your contribution. Left one comment. Also please ensure to sign the DCO

@@ -328,8 +326,7 @@ def __init__(
self._local_scheduling = NodeAffinitySchedulingStrategy(
ray.get_runtime_context().get_node_id(), soft=False
)
Copy link
Contributor

@goutamvenkat-anyscale goutamvenkat-anyscale Jan 12, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@lee1258561 We can't remove this attribute cause there are other dependencies that need it.

What I recommend instead is to call ray.put() on the paths and then have a property based function called _source_paths. That way only the ObjectRef is serialized and not all the paths.

You can look how we do this for _paths in file_based_datasource

Link:

def _paths(self) -> List[str]:
return ray.get(self._paths_ref)

Copy link
Contributor Author

@lee1258561 lee1258561 Feb 3, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I changed the PR based on your suggestion. However, I did not see any other place in this repo referencing _source_paths except the three places that is added in this PR #55978.

Please help me double check if any other places I need to be aware of. Thanks!

@goutamvenkat-anyscale goutamvenkat-anyscale added the @author-action-required The PR author is responsible for the next step. Remove tag to send back to the reviewer. label Jan 13, 2026
@github-actions
Copy link

This pull request has been automatically marked as stale because it has not had
any activity for 14 days. It will be closed in another 14 days if no further activity occurs.
Thank you for your contributions.

You can always ask for help on our discussion forum or Ray's public slack channel.

If you'd like to keep this open, just leave any comment, and the stale label will be removed.

@github-actions github-actions bot added the stale The issue is stale. It will be closed within 7 days unless there are further conversation label Jan 27, 2026
@lee1258561 lee1258561 changed the title Revert "[Data] - Add _source_paths to filedatasources (#57574)" [Data] Store _source_paths in object store rather than self to prevent excessive spilling during read task serialization Feb 3, 2026
@lee1258561 lee1258561 force-pushed the jiunyulee/revert branch 2 times, most recently from 15478ac to 02f68f9 Compare February 3, 2026 08:29
@lee1258561 lee1258561 removed the stale The issue is stale. It will be closed within 7 days unless there are further conversation label Feb 3, 2026
)"

This reverts commit 460bce4.

Signed-off-by: Raymond Lee <lee1258561@gmail.com>
…t ref in self

Signed-off-by: Raymond Lee <lee1258561@gmail.com>
@lee1258561
Copy link
Contributor Author

done signing commit

Signed-off-by: lee1258561 <lee1258561@gmail.com>
@lee1258561 lee1258561 removed the @author-action-required The PR author is responsible for the next step. Remove tag to send back to the reviewer. label Feb 3, 2026
Copy link
Contributor

@goutamvenkat-anyscale goutamvenkat-anyscale left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

lgtm! Thanks for the change

@bveeramani bveeramani merged commit 039dffa into ray-project:master Feb 3, 2026
6 checks passed
rayhhome pushed a commit to rayhhome/ray that referenced this pull request Feb 4, 2026
…t excessive spilling during read task serialization (ray-project#59999)

## Description
This PR reintroducing the issue that A previous PR trying to solve:
ray-project#55978

Specifically, this add a full list of paths to `self` and for Datasource
self is captured every read_task_fn during serialization and causing
this data being duplicated and cause excessive spilling.

We face the similar warning and have spilling behavior:
```
The serialized size of your read function named 'read_task_fn' is 9.7MB. This size relatively large. As a result, Ray might excessively spill objects during execution. To fix this issue, avoid accessing `self` or other large objects in 'read_task_fn'.
```

Revert can solve the issue but this attribute seems to have other
dependancy.

This PR move the _source_paths to object store during init time and only
keep the object ref in self. and introduce a property function to
materialize raw path data when needed

## Related issues
ray-project#55978

## Additional information
> Optional: Add implementation details, API changes, usage examples,
screenshots, etc.

---------

Signed-off-by: Raymond Lee <lee1258561@gmail.com>
Signed-off-by: lee1258561 <lee1258561@gmail.com>
Signed-off-by: Sirui Huang <ray.huang@anyscale.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

community-contribution Contributed by the community data Ray Data-related issues go add ONLY when ready to merge, run all tests

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants